1   package org.apache.lucene.search;
2   
3   /*
4    * Licensed to the Apache Software Foundation (ASF) under one or more
5    * contributor license agreements.  See the NOTICE file distributed with
6    * this work for additional information regarding copyright ownership.
7    * The ASF licenses this file to You under the Apache License, Version 2.0
8    * (the "License"); you may not use this file except in compliance with
9    * the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collection;
23  import java.util.Collections;
24  import java.util.ConcurrentModificationException;
25  import java.util.IdentityHashMap;
26  import java.util.Iterator;
27  import java.util.LinkedHashMap;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Set;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  
33  import org.apache.lucene.index.LeafReader.CoreClosedListener;
34  import org.apache.lucene.index.LeafReaderContext;
35  import org.apache.lucene.index.ReaderUtil;
36  import org.apache.lucene.index.Term;
37  import org.apache.lucene.util.Accountable;
38  import org.apache.lucene.util.Accountables;
39  import org.apache.lucene.util.RamUsageEstimator;
40  import org.apache.lucene.util.RoaringDocIdSet;
41  
42  /**
43   * A {@link QueryCache} that evicts queries using a LRU (least-recently-used)
44   * eviction policy in order to remain under a given maximum size and number of
45   * bytes used.
46   *
47   * This class is thread-safe.
48   *
49   * Note that query eviction runs in linear time with the total number of
50   * segments that have cache entries so this cache works best with
51   * {@link QueryCachingPolicy caching policies} that only cache on "large"
52   * segments, and it is advised to not share this cache across too many indices.
53   *
54   * Typical usage looks like this:
55   * <pre class="prettyprint">
56   *   final int maxNumberOfCachedQueries = 256;
57   *   final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
58   *   // these cache and policy instances can be shared across several queries and readers
59   *   // it is fine to eg. store them into static variables
60   *   final QueryCache queryCache = new LRUQueryCache(maxNumberOfCachedQueries, maxRamBytesUsed);
61   *   final QueryCachingPolicy defaultCachingPolicy = new UsageTrackingQueryCachingPolicy();
62   *
63   *   // ...
64   *
65   *   // Then at search time
66   *   Query myQuery = ...;
67   *   Query myCacheQuery = queryCache.doCache(myQuery, defaultCachingPolicy);
68   *   // myCacheQuery is now a wrapper around the original query that will interact with the cache
69   *   IndexSearcher searcher = ...;
70   *   TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheQuery), 10);
71   * </pre>
72   *
73   * This cache exposes some global statistics ({@link #getHitCount() hit count},
74   * {@link #getMissCount() miss count}, {@link #getCacheSize() number of cache
75   * entries}, {@link #getCacheCount() total number of DocIdSets that have ever
76   * been cached}, {@link #getEvictionCount() number of evicted entries}). In
77   * case you would like to have more fine-grained statistics, such as per-index
78   * or per-query-class statistics, it is possible to override various callbacks:
79   * {@link #onHit}, {@link #onMiss},
80   * {@link #onQueryCache}, {@link #onQueryEviction},
81   * {@link #onDocIdSetCache}, {@link #onDocIdSetEviction} and {@link #onClear}.
82   * It is better to not perform heavy computations in these methods though since
83   * they are called synchronously and under a lock.
84   *
85   * @see QueryCachingPolicy
86   * @lucene.experimental
87   */
88  public class LRUQueryCache implements QueryCache, Accountable {
89  
90    // memory usage of a simple term query
91    static final long QUERY_DEFAULT_RAM_BYTES_USED = 192;
92  
93    static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
94        2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
95        * 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
96  
97    static final long LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY =
98        HASHTABLE_RAM_BYTES_PER_ENTRY
99        + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // previous & next references
100 
101   private final int maxSize;
102   private final long maxRamBytesUsed;
103   // maps queries that are contained in the cache to a singleton so that this
104   // cache does not store several copies of the same query
105   private final Map<Query, Query> uniqueQueries;
106   // The contract between this set and the per-leaf caches is that per-leaf caches
107   // are only allowed to store sub-sets of the queries that are contained in
108   // mostRecentlyUsedQueries. This is why write operations are performed under a lock
109   private final Set<Query> mostRecentlyUsedQueries;
110   private final Map<Object, LeafCache> cache;
111 
112   // these variables are volatile so that we do not need to sync reads
113   // but increments need to be performed under the lock
114   private volatile long ramBytesUsed;
115   private volatile long hitCount;
116   private volatile long missCount;
117   private volatile long cacheCount;
118   private volatile long cacheSize;
119 
120   /**
121    * Create a new instance that will cache at most <code>maxSize</code> queries
122    * with at most <code>maxRamBytesUsed</code> bytes of memory.
123    */
124   public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
125     this.maxSize = maxSize;
126     this.maxRamBytesUsed = maxRamBytesUsed;
127     uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
128     mostRecentlyUsedQueries = uniqueQueries.keySet();
129     cache = new IdentityHashMap<>();
130     ramBytesUsed = 0;
131   }
132 
133   /**
134    * Expert: callback when there is a cache hit on a given query.
135    * Implementing this method is typically useful in order to compute more
136    * fine-grained statistics about the query cache.
137    * @see #onMiss
138    * @lucene.experimental
139    */
140   protected void onHit(Object readerCoreKey, Query query) {
141     hitCount += 1;
142   }
143 
144   /**
145    * Expert: callback when there is a cache miss on a given query.
146    * @see #onHit
147    * @lucene.experimental
148    */
149   protected void onMiss(Object readerCoreKey, Query query) {
150     assert query != null;
151     missCount += 1;
152   }
153 
154   /**
155    * Expert: callback when a query is added to this cache.
156    * Implementing this method is typically useful in order to compute more
157    * fine-grained statistics about the query cache.
158    * @see #onQueryEviction
159    * @lucene.experimental
160    */
161   protected void onQueryCache(Query query, long ramBytesUsed) {
162     this.ramBytesUsed += ramBytesUsed;
163   }
164 
165   /**
166    * Expert: callback when a query is evicted from this cache.
167    * @see #onQueryCache
168    * @lucene.experimental
169    */
170   protected void onQueryEviction(Query query, long ramBytesUsed) {
171     this.ramBytesUsed -= ramBytesUsed;
172   }
173 
174   /**
175    * Expert: callback when a {@link DocIdSet} is added to this cache.
176    * Implementing this method is typically useful in order to compute more
177    * fine-grained statistics about the query cache.
178    * @see #onDocIdSetEviction
179    * @lucene.experimental
180    */
181   protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
182     cacheSize += 1;
183     cacheCount += 1;
184     this.ramBytesUsed += ramBytesUsed;
185   }
186 
187   /**
188    * Expert: callback when one or more {@link DocIdSet}s are removed from this
189    * cache.
190    * @see #onDocIdSetCache
191    * @lucene.experimental
192    */
193   protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
194     this.ramBytesUsed -= sumRamBytesUsed;
195     cacheSize -= numEntries;
196   }
197 
198   /**
199    * Expert: callback when the cache is completely cleared.
200    * @lucene.experimental
201    */
202   protected void onClear() {
203     ramBytesUsed = 0;
204     cacheSize = 0;
205   }
206 
207   /** Whether evictions are required. */
208   boolean requiresEviction() {
209     final int size = mostRecentlyUsedQueries.size();
210     if (size == 0) {
211       return false;
212     } else {
213       return size > maxSize || ramBytesUsed() > maxRamBytesUsed;
214     }
215   }
216 
217   synchronized DocIdSet get(Query key, LeafReaderContext context) {
218     assert key.getBoost() == 1f;
219     assert key instanceof BoostQuery == false;
220     assert key instanceof ConstantScoreQuery == false;
221     final Object readerKey = context.reader().getCoreCacheKey();
222     final LeafCache leafCache = cache.get(readerKey);
223     if (leafCache == null) {
224       onMiss(readerKey, key);
225       return null;
226     }
227     // this get call moves the query to the most-recently-used position
228     final Query singleton = uniqueQueries.get(key);
229     if (singleton == null) {
230       onMiss(readerKey, key);
231       return null;
232     }
233     final DocIdSet cached = leafCache.get(singleton);
234     if (cached == null) {
235       onMiss(readerKey, singleton);
236     } else {
237       onHit(readerKey, singleton);
238     }
239     return cached;
240   }
241 
242   synchronized void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
243     // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
244     // we don't want to have user-provided queries as keys in our cache since queries are mutable
245     assert query instanceof BoostQuery == false;
246     assert query instanceof ConstantScoreQuery == false;
247     assert query.getBoost() == 1f;
248     Query singleton = uniqueQueries.get(query);
249     if (singleton == null) {
250       uniqueQueries.put(query, query);
251       onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query));
252     } else {
253       query = singleton;
254     }
255     final Object key = context.reader().getCoreCacheKey();
256     LeafCache leafCache = cache.get(key);
257     if (leafCache == null) {
258       leafCache = new LeafCache(key);
259       final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
260       ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
261       assert previous == null;
262       // we just created a new leaf cache, need to register a close listener
263       context.reader().addCoreClosedListener(new CoreClosedListener() {
264         @Override
265         public void onClose(Object ownerCoreCacheKey) {
266           clearCoreCacheKey(ownerCoreCacheKey);
267         }
268       });
269     }
270     leafCache.putIfAbsent(query, set);
271     evictIfNecessary();
272   }
273 
274   synchronized void evictIfNecessary() {
275     // under a lock to make sure that mostRecentlyUsedQueries and cache keep sync'ed
276     if (requiresEviction()) {
277       
278       Iterator<Query> iterator = mostRecentlyUsedQueries.iterator();
279       do {
280         final Query query = iterator.next();
281         final int size = mostRecentlyUsedQueries.size();
282         iterator.remove();
283         if (size == mostRecentlyUsedQueries.size()) {
284           // size did not decrease, because the hash of the query changed since it has been
285           // put into the cache
286           throw new ConcurrentModificationException("Removal from the cache failed! This " +
287               "is probably due to a query which has been modified after having been put into " +
288               " the cache or a badly implemented clone(). Query class: [" + query.getClass() +
289               "], query: [" + query + "]");
290         }
291         onEviction(query);
292       } while (iterator.hasNext() && requiresEviction());
293     }
294   }
295 
296   /**
297    * Remove all cache entries for the given core cache key.
298    */
299   public synchronized void clearCoreCacheKey(Object coreKey) {
300     final LeafCache leafCache = cache.remove(coreKey);
301     if (leafCache != null) {
302       ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
303       onDocIdSetEviction(coreKey, leafCache.cache.size(), leafCache.ramBytesUsed);
304     }
305   }
306 
307   /**
308    * Remove all cache entries for the given query.
309    */
310   public synchronized void clearQuery(Query query) {
311     final Query singleton = uniqueQueries.remove(query);
312     if (singleton != null) {
313       onEviction(singleton);
314     }
315   }
316 
317   private void onEviction(Query singleton) {
318     onQueryEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
319     for (LeafCache leafCache : cache.values()) {
320       leafCache.remove(singleton);
321     }
322   }
323 
324   /**
325    * Clear the content of this cache.
326    */
327   public synchronized void clear() {
328     cache.clear();
329     mostRecentlyUsedQueries.clear();
330     onClear();
331   }
332 
333   // pkg-private for testing
334   synchronized void assertConsistent() {
335     if (requiresEviction()) {
336       throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size()
337           + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
338     }
339     for (LeafCache leafCache : cache.values()) {
340       Set<Query> keys = Collections.newSetFromMap(new IdentityHashMap<Query, Boolean>());
341       keys.addAll(leafCache.cache.keySet());
342       keys.removeAll(mostRecentlyUsedQueries);
343       if (!keys.isEmpty()) {
344         throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
345       }
346     }
347     long recomputedRamBytesUsed =
348           HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
349         + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size();
350     for (Query query : mostRecentlyUsedQueries) {
351       recomputedRamBytesUsed += ramBytesUsed(query);
352     }
353     for (LeafCache leafCache : cache.values()) {
354       recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
355       for (DocIdSet set : leafCache.cache.values()) {
356         recomputedRamBytesUsed += set.ramBytesUsed();
357       }
358     }
359     if (recomputedRamBytesUsed != ramBytesUsed) {
360       throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
361     }
362 
363     long recomputedCacheSize = 0;
364     for (LeafCache leafCache : cache.values()) {
365       recomputedCacheSize += leafCache.cache.size();
366     }
367     if (recomputedCacheSize != getCacheSize()) {
368       throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize);
369     }
370   }
371 
372   // pkg-private for testing
373   // return the list of cached queries in LRU order
374   synchronized List<Query> cachedQueries() {
375     return new ArrayList<>(mostRecentlyUsedQueries);
376   }
377 
378   @Override
379   public Weight doCache(Weight weight, QueryCachingPolicy policy) {
380     while (weight instanceof CachingWrapperWeight) {
381       weight = ((CachingWrapperWeight) weight).in;
382     }
383 
384     return new CachingWrapperWeight(weight, policy);
385   }
386 
387   @Override
388   public long ramBytesUsed() {
389     return ramBytesUsed;
390   }
391 
392   @Override
393   public Collection<Accountable> getChildResources() {
394     synchronized (this) {
395       return Accountables.namedAccountables("segment", cache);
396     }
397   }
398 
399   /**
400    * Return the number of bytes used by the given query. The default
401    * implementation returns {@link Accountable#ramBytesUsed()} if the query
402    * implements {@link Accountable} and <code>1024</code> otherwise.
403    */
404   protected long ramBytesUsed(Query query) {
405     if (query instanceof Accountable) {
406       return ((Accountable) query).ramBytesUsed();
407     }
408     return QUERY_DEFAULT_RAM_BYTES_USED;
409   }
410 
411   /**
412    * Default cache implementation: uses {@link RoaringDocIdSet}.
413    */
414   protected DocIdSet cacheImpl(BulkScorer scorer, int maxDoc) throws IOException {
415     final RoaringDocIdSet.Builder builder = new RoaringDocIdSet.Builder(maxDoc);
416     scorer.score(new LeafCollector() {
417 
418       @Override
419       public void setScorer(Scorer scorer) throws IOException {}
420 
421       @Override
422       public void collect(int doc) throws IOException {
423         builder.add(doc);
424       }
425 
426     }, null);
427     return builder.build();
428   }
429 
430   /**
431    * Return the total number of times that a {@link Query} has been looked up
432    * in this {@link QueryCache}. Note that this number is incremented once per
433    * segment so running a cached query only once will increment this counter
434    * by the number of segments that are wrapped by the searcher.
435    * Note that by definition, {@link #getTotalCount()} is the sum of
436    * {@link #getHitCount()} and {@link #getMissCount()}.
437    * @see #getHitCount()
438    * @see #getMissCount()
439    */
440   public final long getTotalCount() {
441     return getHitCount() + getMissCount();
442   }
443 
444   /**
445    * Over the {@link #getTotalCount() total} number of times that a query has
446    * been looked up, return how many times a cached {@link DocIdSet} has been
447    * found and returned.
448    * @see #getTotalCount()
449    * @see #getMissCount()
450    */
451   public final long getHitCount() {
452     return hitCount;
453   }
454 
455   /**
456    * Over the {@link #getTotalCount() total} number of times that a query has
457    * been looked up, return how many times this query was not contained in the
458    * cache.
459    * @see #getTotalCount()
460    * @see #getHitCount()
461    */
462   public final long getMissCount() {
463     return missCount;
464   }
465 
466   /**
467    * Return the total number of {@link DocIdSet}s which are currently stored
468    * in the cache.
469    * @see #getCacheCount()
470    * @see #getEvictionCount()
471    */
472   public final long getCacheSize() {
473     return cacheSize;
474   }
475 
476   /**
477    * Return the total number of cache entries that have been generated and put
478    * in the cache. It is highly desirable to have a {@link #getHitCount() hit
479    * count} that is much higher than the {@link #getCacheCount() cache count}
480    * as the opposite would indicate that the query cache makes efforts in order
481    * to cache queries but then they do not get reused.
482    * @see #getCacheSize()
483    * @see #getEvictionCount()
484    */
485   public final long getCacheCount() {
486     return cacheCount;
487   }
488 
489   /**
490    * Return the number of cache entries that have been removed from the cache
491    * either in order to stay under the maximum configured size/ram usage, or
492    * because a segment has been closed. High numbers of evictions might mean
493    * that queries are not reused or that the {@link QueryCachingPolicy
494    * caching policy} caches too aggressively on NRT segments which get merged
495    * early.
496    * @see #getCacheCount()
497    * @see #getCacheSize()
498    */
499   public final long getEvictionCount() {
500     return getCacheCount() - getCacheSize();
501   }
502 
503   // this class is not thread-safe, everything but ramBytesUsed needs to be called under a lock
504   private class LeafCache implements Accountable {
505 
506     private final Object key;
507     private final Map<Query, DocIdSet> cache;
508     private volatile long ramBytesUsed;
509 
510     LeafCache(Object key) {
511       this.key = key;
512       cache = new IdentityHashMap<>();
513       ramBytesUsed = 0;
514     }
515 
516     private void onDocIdSetCache(long ramBytesUsed) {
517       this.ramBytesUsed += ramBytesUsed;
518       LRUQueryCache.this.onDocIdSetCache(key, ramBytesUsed);
519     }
520 
521     private void onDocIdSetEviction(long ramBytesUsed) {
522       this.ramBytesUsed -= ramBytesUsed;
523       LRUQueryCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
524     }
525 
526     DocIdSet get(Query query) {
527       assert query instanceof BoostQuery == false;
528       assert query instanceof ConstantScoreQuery == false;
529       assert query.getBoost() == 1f;
530       return cache.get(query);
531     }
532 
533     void putIfAbsent(Query query, DocIdSet set) {
534       assert query instanceof BoostQuery == false;
535       assert query instanceof ConstantScoreQuery == false;
536       assert query.getBoost() == 1f;
537       if (cache.containsKey(query) == false) {
538         cache.put(query, set);
539         // the set was actually put
540         onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
541       }
542     }
543 
544     void remove(Query query) {
545       assert query instanceof BoostQuery == false;
546       assert query instanceof ConstantScoreQuery == false;
547       assert query.getBoost() == 1f;
548       DocIdSet removed = cache.remove(query);
549       if (removed != null) {
550         onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed());
551       }
552     }
553 
554     @Override
555     public long ramBytesUsed() {
556       return ramBytesUsed;
557     }
558 
559     @Override
560     public Collection<Accountable> getChildResources() {
561       return Collections.emptyList();
562     }
563   }
564 
565   private class CachingWrapperWeight extends ConstantScoreWeight {
566 
567     private final Weight in;
568     private final QueryCachingPolicy policy;
569     // we use an AtomicBoolean because Weight.scorer may be called from multiple
570     // threads when IndexSearcher is created with threads
571     private final AtomicBoolean used;
572 
573     CachingWrapperWeight(Weight in, QueryCachingPolicy policy) {
574       super(in.getQuery());
575       this.in = in;
576       this.policy = policy;
577       used = new AtomicBoolean(false);
578     }
579 
580     @Override
581     public void extractTerms(Set<Term> terms) {
582       in.extractTerms(terms);
583     }
584 
585     private boolean cacheEntryHasReasonableWorstCaseSize(int maxDoc) {
586       // The worst-case (dense) is a bit set which needs one bit per document
587       final long worstCaseRamUsage = maxDoc / 8;
588       final long totalRamAvailable = maxRamBytesUsed;
589       // Imagine the worst-case that a cache entry is large than the size of
590       // the cache: not only will this entry be trashed immediately but it
591       // will also evict all current entries from the cache. For this reason
592       // we only cache on an IndexReader if we have available room for
593       // 5 different filters on this reader to avoid excessive trashing
594       return worstCaseRamUsage * 5 < totalRamAvailable;
595     }
596 
597     private DocIdSet cache(LeafReaderContext context) throws IOException {
598       final BulkScorer scorer = in.bulkScorer(context);
599       if (scorer == null) {
600         return DocIdSet.EMPTY;
601       } else {
602         return cacheImpl(scorer, context.reader().maxDoc());
603       }
604     }
605 
606     private boolean shouldCache(LeafReaderContext context) throws IOException {
607       return cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
608           && policy.shouldCache(in.getQuery(), context);
609     }
610 
611     @Override
612     public Scorer scorer(LeafReaderContext context) throws IOException {
613       if (used.compareAndSet(false, true)) {
614         policy.onUse(getQuery());
615       }
616       DocIdSet docIdSet = get(in.getQuery(), context);
617       if (docIdSet == null) {
618         if (shouldCache(context)) {
619           docIdSet = cache(context);
620           putIfAbsent(in.getQuery(), context, docIdSet);
621         } else {
622           return in.scorer(context);
623         }
624       }
625 
626       assert docIdSet != null;
627       if (docIdSet == DocIdSet.EMPTY) {
628         return null;
629       }
630       final DocIdSetIterator disi = docIdSet.iterator();
631       if (disi == null) {
632         return null;
633       }
634 
635       return new ConstantScoreScorer(this, 0f, disi);
636     }
637 
638     @Override
639     public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
640       if (used.compareAndSet(false, true)) {
641         policy.onUse(getQuery());
642       }
643       DocIdSet docIdSet = get(in.getQuery(), context);
644       if (docIdSet == null) {
645         if (shouldCache(context)) {
646           docIdSet = cache(context);
647           putIfAbsent(in.getQuery(), context, docIdSet);
648         } else {
649           return in.bulkScorer(context);
650         }
651       }
652 
653       assert docIdSet != null;
654       if (docIdSet == DocIdSet.EMPTY) {
655         return null;
656       }
657       final DocIdSetIterator disi = docIdSet.iterator();
658       if (disi == null) {
659         return null;
660       }
661 
662       return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, disi));
663     }
664 
665   }
666 }